92707c
@@ -12230,7 +12230,7 @@
private boolean canHandleQuery(QB qbToChk, boolean topLevelQB) {
     private RelOptSchema                                  relOptSchema;
     private SemanticException                             semanticException;
     private Map<String, PrunedPartitionList>              partitionCache;
-    private AtomicInteger                                 noColsMissingStats = new AtomicInteger(0);
+    private final AtomicInteger                                 noColsMissingStats = new AtomicInteger(0);
     List<FieldSchema>                                     topLevelFieldSchema;
 
     // TODO: Do we need to keep track of RR, ColNameToPosMap for every op or
@@ -12808,6 +12808,14 @@
private RelNode genTableLogicalPlan(String tableAlias, QB qb) throws SemanticExc
 
     private RelNode genFilterRelNode(ASTNode filterExpr, RelNode srcRel) throws SemanticException {
       ExprNodeDesc filterCondn = genExprNodeDesc(filterExpr, relToHiveRR.get(srcRel));
+      if (filterCondn instanceof ExprNodeConstantDesc &&
+        !filterCondn.getTypeString().equals(serdeConstants.BOOLEAN_TYPE_NAME)) {
+        // queries like select * from t1 where 'foo';
+        // Optiq's rule PushFilterThroughProject chokes on it. Arguably, we can insert a cast to
+        // boolean in such cases, but since Postgres, Oracle and MS SQL server fail on compile time
+        // for such queries, its an arcane corner case, not worth of adding that complexity.
+        throw new OptiqSemanticException("Filter expression with non-boolean return type.");
+      }
       ImmutableMap<String, Integer> hiveColNameOptiqPosMap = this.relToHiveColNameOptiqPosMap
           .get(srcRel);
       RexNode convertedFilterExpr = new RexNodeConverter(cluster, srcRel.getRowType(),
